9a297e
@@ -128,14 +128,15 @@
public class HTable implements HTableInterface {
     if (maxThreads == 0) {
       maxThreads = 1; // is there a better default?
     }
+    int corePoolSize = conf.getInt("hbase.htable.threads.coresize", 1);
     long keepAliveTime = conf.getLong("hbase.htable.threads.keepalivetime", 60);
 
     // Using the "direct handoff" approach, new threads will only be created
     // if it is necessary and will grow unbounded. This could be bad but in HCM
     // we only create as many Runnables as there are region servers. It means
     // it also scales when new region servers are added.
-    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, TimeUnit.SECONDS,
-        new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
+    ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime,
+      TimeUnit.SECONDS, new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
     pool.allowCoreThreadTimeOut(true);
     return pool;
   }
